fc2464
@@ -47,6 +47,21 @@
public abstract class AbstractDataStreamWriter extends OutputStoreObjectSupport
 
 	private int maxOpenAttempts = DEFAULT_MAX_OPEN_ATTEMPTS;
 
+	/**
+	 * We use this jvm level lock in this class to guard against one
+	 * scenario. When we try to create a stream a check is first done
+	 * if path exists and then we create a stream. This sync lock would
+	 * not make sense on a global level because exist()/create() is never
+	 * atomic but we want to do this within a jvm. Some distros have a little
+	 * different functionality in cases when same leaseholder is trying to
+	 * re-create a stream with already open file. i.e. in cdh this operation
+	 * on default takes 5 minutes while vanilla hadoop fails immediately.
+	 * We minimise this risk within a jvm in a way that same leaseholder
+	 * would not try to use same path to create a stream. In different jvm's
+	 * error handling is different because of different leaseholders.
+	 */
+	private final static Object lock = new Object();
+
 	/**
 	 * Instantiates a new abstract data stream writer.
 	 *
@@ -89,16 +104,30 @@
public abstract class AbstractDataStreamWriter extends OutputStoreObjectSupport
 		Path p = null;
 		FSDataOutputStream wout = null;
 		int openAttempt = 0;
+
 		do {
+			boolean rollStrategies = false;
 			try {
 				p = getResolvedPath();
-				if (isAppendable() && p.getFileSystem(getConfiguration()).exists(p)) {
-					wout = fs.append(p);
-				} else {
-					wout = fs.create(p, isOverwrite());
+				synchronized (lock) {
+					boolean exists = fs.exists(p);
+					if (isAppendable() && exists) {
+						wout = fs.append(p);
+						break;
+					} else if (!isOverwrite() && exists) {
+						// don't rely on error to roll.
+						// check notes for lock object.
+						rollStrategies = true;
+					} else {
+						wout = fs.create(p, isOverwrite());
+						break;
+					}
 				}
-				break;
 			} catch (Exception e) {
+				rollStrategies = true;
+			}
+
+			if (rollStrategies) {
 				getOutputContext().rollStrategies();
 			}
 
